8a06b9
@@ -58,6 +58,7 @@
 import javax.jdo.JDOException;
 
 import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -2937,8 +2938,8 @@
public Partition exchange_partition(Map<String, String> partitionSpecs,
           Warehouse.makePartName(partitionKeysPresent, partValsPresent));
       Path destPath = new Path(destinationTable.getSd().getLocation(),
           Warehouse.makePartName(partitionKeysPresent, partValsPresent));
+      List<Partition> destPartitions = new ArrayList<Partition>();
       try {
-        List<Partition> destPartitions = new ArrayList<Partition>();
         for (Partition partition: partitionsToExchange) {
           Partition destPartition = new Partition(partition);
           destPartition.setDbName(destDbName);
@@ -2962,6 +2963,12 @@
public Partition exchange_partition(Map<String, String> partitionSpecs,
          * once https://issues.apache.org/jira/browse/HDFS-3370 is done
          */
         pathCreated = wh.renameDir(sourcePath, destPath);
+
+        // Setting success to false to make sure that if the listener fails, rollback happens.
+        success = false;
+        fireMetaStoreExchangePartitionEvent(sourceTable, partitionsToExchange,
+            destinationTable, destPartitions, transactionalListeners, true);
+
         success = ms.commitTransaction();
         return destPartitions;
       } finally {
@@ -2970,6 +2977,35 @@
public Partition exchange_partition(Map<String, String> partitionSpecs,
           if (pathCreated) {
             wh.renameDir(destPath, sourcePath);
           }
+
+          fireMetaStoreExchangePartitionEvent(sourceTable, partitionsToExchange,
+              destinationTable, destPartitions, listeners, success);
+        }
+      }
+    }
+
+    private void fireMetaStoreExchangePartitionEvent(Table sourceTable,
+        List<Partition> partitionsToExchange, Table destinationTable,
+        List<Partition> destPartitions,
+        List<MetaStoreEventListener> eventListeners,
+        boolean status) throws MetaException {
+      if (sourceTable != null && destinationTable != null
+          && !CollectionUtils.isEmpty(partitionsToExchange)
+          && !CollectionUtils.isEmpty(destPartitions)) {
+        if (eventListeners.size() > 0) {
+          AddPartitionEvent addPartitionEvent =
+              new AddPartitionEvent(destinationTable, destPartitions, status, this);
+          for (MetaStoreEventListener eventListener : eventListeners) {
+            eventListener.onAddPartition(addPartitionEvent);
+          }
+
+          for (Partition partition : partitionsToExchange) {
+            DropPartitionEvent dropPartitionEvent =
+                new DropPartitionEvent(sourceTable, partition, true, status, this);
+            for (MetaStoreEventListener eventListener : eventListeners) {
+              eventListener.onDropPartition(dropPartitionEvent);
+            }
+          }
         }
       }
     }
